Skip to content

[fix][broker] Fix compacted read could be stuck forever or message loss due to cursor mark delete#25998

Merged
BewareMyPower merged 6 commits into
apache:masterfrom
BewareMyPower:bewaremypower/fix-read-compacted-topic-stuck
Jun 12, 2026
Merged

[fix][broker] Fix compacted read could be stuck forever or message loss due to cursor mark delete#25998
BewareMyPower merged 6 commits into
apache:masterfrom
BewareMyPower:bewaremypower/fix-read-compacted-topic-stuck

Conversation

@BewareMyPower

@BewareMyPower BewareMyPower commented Jun 11, 2026

Copy link
Copy Markdown
Contributor

Motivation

We observed the system topic reader encountered a typical bug case: hasMessageAvailable() returns true but readNext() is not able to read any message. It causes all topics in the same namespace fail to load.

After digging into details, I've figured out the cause for this incident. During the incident, there are two unload operations on bundles of the same namespace in short time:

  1. The 1st bundle unloaded a user topic, then the topic loading triggered the read on __change_events topic
  2. The system topic reader started to read from earliest, then it read a message from broker-0, the message id is A:2
  3. The 2nd bundle unloaded the __change_events topic from broker-0 to broker-1
  4. The system topic reader then reconnected to broker-1, with A:2 as the start message id.

Then the reader was stuck. From the heap dump, I found the following important info from the dispatcher:

isFirstRead: true
activeConsumer:
  msgOutCounter: 0
  messagePermits: 1000
  startMessageId: "A:2"
  msgOutCounter: 0
  messageAckCounter: 1
cursor:
  ledger:
    ledgers:
    - {"ledgerId":B,"entries":0}
    lastConfirmedEntry: "B:-1"
  entriesReadCount: 0
  readPosition: "B:0"
  waitingReadOp:
    entries: []
    readPosition: "B:0"
    nextReadPosition: "B:0"

As you can see, all entries are compacted (the original ledger id is A), the managed ledger was just empty with an empty ledger whose id is B.

From the logs, we can see the cursor's initial mark delete position and read position is different:

ackPos=A:1, readPos=A:2

But the ackPos is A:2 in the heap dump, and readPos is B:0. I also checked the topic compaction context's cache and dumped the compacted ledger to verify there is even no findStartPoint call on the compacted ledger.

The root cause is:

  1. When the reader received a message from broker-0, it stored the message ID A:2 to PersistentAcknowledgmentsGroupingTracker#lastCumulativeAck via acknowledgeCumulativeAsync.
  2. After 100ms (the default acknowledgmentGroupTime), the reader had already switched the connection to broker-1, then it flushed the ACK request to broker-1
  3. broker-1 received the ACK request and simply called cursor.asyncMarkDelete(A:2).

cursor.asyncMarkDelete(position, mergeCursorProperties(properties),

The mark delete position was simply set to A:2:

However, the read position was advanced to the next valid position (B:0) in the managed ledger:

Position newReadPosition = ledger.getNextValidPosition(markDeletePosition);

while the compaction horizon is just B:-1, which is behind the new read position:

|| readPosition.compareTo(
lastCompactedPosition.getLedgerId(), lastCompactedPosition.getEntryId()) > 0) {

and the read operation went through the managed cursor directly and would be stuck forever.

Modifications

When a subscription has a compacted consumer, if the acknowledged position's ledger id does not exist in the managed ledger, just do nothing. Otherwise, even if the next valid ledger has messages to read, the compacted entries could be skipped, which leads to message loss.

Verifying this change

testReceiveAckAfterReconnectionOnEmptyLedger covers the case exactly.

Others

Consumer does not have this issue, even if it's non-durable, because consumer's startMessageId is null, so in asyncReadCompactedEntries, in the first read, readFromEarliest is always true.

@BewareMyPower BewareMyPower self-assigned this Jun 11, 2026
@BewareMyPower BewareMyPower changed the title [fix][broker] Fix compacted read could be stuck forever or message loss due to delayed acknowledgment [fix][broker] Fix compacted read could be stuck forever or message loss due to cursor mark delete Jun 11, 2026
@codelipenghui codelipenghui added this to the 5.0.0-M1 milestone Jun 11, 2026
@BewareMyPower

Copy link
Copy Markdown
Contributor Author

PublishRateLimiterOverconsumingTest looks like a flaky tes

@BewareMyPower BewareMyPower marked this pull request as draft June 11, 2026 11:33
@BewareMyPower

Copy link
Copy Markdown
Contributor Author

After removing conf.setSystemTopicEnabled(false), the test will be hard to reproduce the issue, I will improve the test

@BewareMyPower

Copy link
Copy Markdown
Contributor Author

Oh I've figured out the reason, it's because both system topic reader and the normal reader have reader in the subscription name

@BewareMyPower BewareMyPower marked this pull request as ready for review June 11, 2026 12:19

@void-ptr974 void-ptr974 left a comment

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the fix! The updated change looks good to me.

@BewareMyPower BewareMyPower merged commit 7a9fefb into apache:master Jun 12, 2026
81 of 83 checks passed
@BewareMyPower BewareMyPower deleted the bewaremypower/fix-read-compacted-topic-stuck branch June 12, 2026 03:34
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants